1   package org.apache.lucene.search;
2   
3   /*
4    * Licensed to the Apache Software Foundation (ASF) under one or more
5    * contributor license agreements.  See the NOTICE file distributed with
6    * this work for additional information regarding copyright ownership.
7    * The ASF licenses this file to You under the Apache License, Version 2.0
8    * (the "License"); you may not use this file except in compliance with
9    * the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  
20  import java.io.IOException;
21  import java.util.ArrayList;
22  import java.util.Collection;
23  import java.util.Collections;
24  import java.util.IdentityHashMap;
25  import java.util.Iterator;
26  import java.util.LinkedHashMap;
27  import java.util.List;
28  import java.util.Map;
29  import java.util.Set;
30  
31  import org.apache.lucene.index.LeafReader;
32  import org.apache.lucene.index.LeafReader.CoreClosedListener;
33  import org.apache.lucene.index.LeafReaderContext;
34  import org.apache.lucene.util.Accountable;
35  import org.apache.lucene.util.Accountables;
36  import org.apache.lucene.util.Bits;
37  import org.apache.lucene.util.RamUsageEstimator;
38  import org.apache.lucene.util.RoaringDocIdSet;
39  
40  /**
41   * A {@link FilterCache} that evicts filters using a LRU (least-recently-used)
42   * eviction policy in order to remain under a given maximum size and number of
43   * bytes used.
44   *
45   * This class is thread-safe.
46   *
47   * Note that filter eviction runs in linear time with the total number of
48   * segments that have cache entries so this cache works best with
49   * {@link FilterCachingPolicy caching policies} that only cache on "large"
50   * segments, and it is advised to not share this cache across too many indices.
51   *
52   * Typical usage looks like this:
53   * <pre class="prettyprint">
54   *   final int maxNumberOfCachedFilters = 256;
55   *   final long maxRamBytesUsed = 50 * 1024L * 1024L; // 50MB
56   *   // these cache and policy instances can be shared across several filters and readers
57   *   // it is fine to eg. store them into static variables
58   *   final FilterCache filterCache = new LRUFilterCache(maxNumberOfCachedFilters, maxRamBytesUsed);
59   *   final FilterCachingPolicy defaultCachingPolicy = new UsageTrackingFilterCachingPolicy();
60   *   
61   *   // ...
62   *   
63   *   // Then at search time
64   *   Filter myFilter = ...;
65   *   Filter myCacheFilter = filterCache.doCache(myFilter, defaultCachingPolicy);
66   *   // myCacheFilter is now a wrapper around the original filter that will interact with the cache
67   *   IndexSearcher searcher = ...;
68   *   TopDocs topDocs = searcher.search(new ConstantScoreQuery(myCacheFilter), 10);
69   * </pre>
70   *
71   * This cache exposes some global statistics ({@link #getHitCount() hit count},
72   * {@link #getMissCount() miss count}, {@link #getCacheSize() number of cache
73   * entries}, {@link #getCacheCount() total number of DocIdSets that have ever
74   * been cached}, {@link #getEvictionCount() number of evicted entries}). In
75   * case you would like to have more fine-grained statistics, such as per-index
76   * or per-filter-class statistics, it is possible to override various callbacks:
77   * {@link #onHit}, {@link #onMiss},
78   * {@link #onFilterCache}, {@link #onFilterEviction},
79   * {@link #onDocIdSetCache}, {@link #onDocIdSetEviction} and {@link #onClear}.
80   * It is better to not perform heavy computations in these methods though since
81   * they are called synchronously and under a lock.
82   *
83   * @see FilterCachingPolicy
84   * @lucene.experimental
85   * @deprecated Use {@link LRUQueryCache} instead
86   */
87  @Deprecated
88  public class LRUFilterCache implements FilterCache, Accountable {
89  
90    // memory usage of a simple query-wrapper filter around a term query
91    static final long FILTER_DEFAULT_RAM_BYTES_USED = 216;
92  
93    static final long HASHTABLE_RAM_BYTES_PER_ENTRY =
94        2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF // key + value
95        * 2; // hash tables need to be oversized to avoid collisions, assume 2x capacity
96  
97    static final long LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY =
98        HASHTABLE_RAM_BYTES_PER_ENTRY
99        + 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF; // previous & next references
100 
101   private final int maxSize;
102   private final long maxRamBytesUsed;
103   // maps filters that are contained in the cache to a singleton so that this
104   // cache does not store several copies of the same filter
105   private final Map<Filter, Filter> uniqueFilters;
106   // The contract between this set and the per-leaf caches is that per-leaf caches
107   // are only allowed to store sub-sets of the filters that are contained in
108   // mostRecentlyUsedFilters. This is why write operations are performed under a lock
109   private final Set<Filter> mostRecentlyUsedFilters;
110   private final Map<Object, LeafCache> cache;
111 
112   // these variables are volatile so that we do not need to sync reads
113   // but increments need to be performed under the lock
114   private volatile long ramBytesUsed;
115   private volatile long hitCount;
116   private volatile long missCount;
117   private volatile long cacheCount;
118   private volatile long cacheSize;
119 
120   /**
121    * Create a new instance that will cache at most <code>maxSize</code> filters
122    * with at most <code>maxRamBytesUsed</code> bytes of memory.
123    */
124   public LRUFilterCache(int maxSize, long maxRamBytesUsed) {
125     this.maxSize = maxSize;
126     this.maxRamBytesUsed = maxRamBytesUsed;
127     uniqueFilters = new LinkedHashMap<Filter, Filter>(16, 0.75f, true);
128     mostRecentlyUsedFilters = uniqueFilters.keySet();
129     cache = new IdentityHashMap<>();
130     ramBytesUsed = 0;
131   }
132 
133   /**
134    * Expert: callback when there is a cache hit on a given filter.
135    * Implementing this method is typically useful in order to compute more
136    * fine-grained statistics about the filter cache.
137    * @see #onMiss
138    * @lucene.experimental
139    */
140   protected void onHit(Object readerCoreKey, Filter filter) {
141     hitCount += 1;
142   }
143 
144   /**
145    * Expert: callback when there is a cache miss on a given filter.
146    * @see #onHit
147    * @lucene.experimental
148    */
149   protected void onMiss(Object readerCoreKey, Filter filter) {
150     assert filter != null;
151     missCount += 1;
152   }
153 
154   /**
155    * Expert: callback when a filter is added to this cache.
156    * Implementing this method is typically useful in order to compute more
157    * fine-grained statistics about the filter cache.
158    * @see #onFilterEviction
159    * @lucene.experimental
160    */
161   protected void onFilterCache(Filter filter, long ramBytesUsed) {
162     this.ramBytesUsed += ramBytesUsed;
163   }
164 
165   /**
166    * Expert: callback when a filter is evicted from this cache.
167    * @see #onFilterCache
168    * @lucene.experimental
169    */
170   protected void onFilterEviction(Filter filter, long ramBytesUsed) {
171     this.ramBytesUsed -= ramBytesUsed;
172   }
173 
174   /**
175    * Expert: callback when a {@link DocIdSet} is added to this cache.
176    * Implementing this method is typically useful in order to compute more
177    * fine-grained statistics about the filter cache.
178    * @see #onDocIdSetEviction
179    * @lucene.experimental
180    */
181   protected void onDocIdSetCache(Object readerCoreKey, long ramBytesUsed) {
182     cacheSize += 1;
183     cacheCount += 1;
184     this.ramBytesUsed += ramBytesUsed;
185   }
186   
187   /**
188    * Expert: callback when one or more {@link DocIdSet}s are removed from this
189    * cache.
190    * @see #onDocIdSetCache
191    * @lucene.experimental
192    */
193   protected void onDocIdSetEviction(Object readerCoreKey, int numEntries, long sumRamBytesUsed) {
194     this.ramBytesUsed -= sumRamBytesUsed;
195     cacheSize -= numEntries;
196   }
197 
198   /**
199    * Expert: callback when the cache is completely cleared.
200    * @lucene.experimental
201    */
202   protected void onClear() {
203     ramBytesUsed = 0;
204     cacheSize = 0;
205   }
206 
207   /** Whether evictions are required. */
208   boolean requiresEviction() {
209     final int size = mostRecentlyUsedFilters.size();
210     if (size == 0) {
211       return false;
212     } else {
213       return size > maxSize || ramBytesUsed() > maxRamBytesUsed;
214     }
215   }
216 
217   synchronized DocIdSet get(Filter filter, LeafReaderContext context) {
218     final Object readerKey = context.reader().getCoreCacheKey();
219     final LeafCache leafCache = cache.get(readerKey);
220     if (leafCache == null) {
221       onMiss(readerKey, filter);
222       return null;
223     }
224     // this get call moves the filter to the most-recently-used position
225     final Filter singleton = uniqueFilters.get(filter);
226     if (singleton == null) {
227       onMiss(readerKey, filter);
228       return null;
229     }
230     final DocIdSet cached = leafCache.get(singleton);
231     if (cached == null) {
232       onMiss(readerKey, singleton);
233     } else {
234       onHit(readerKey, singleton);
235     }
236     return cached;
237   }
238 
239   synchronized void putIfAbsent(Filter filter, LeafReaderContext context, DocIdSet set) {
240     // under a lock to make sure that mostRecentlyUsedFilters and cache remain sync'ed
241     assert set.isCacheable();
242     Filter singleton = uniqueFilters.get(filter);
243     if (singleton == null) {
244       uniqueFilters.put(filter, filter);
245       onFilterCache(singleton, LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(filter));
246     } else {
247       filter = singleton;
248     }
249     final Object key = context.reader().getCoreCacheKey();
250     LeafCache leafCache = cache.get(key);
251     if (leafCache == null) {
252       leafCache = new LeafCache(key);
253       final LeafCache previous = cache.put(context.reader().getCoreCacheKey(), leafCache);
254       ramBytesUsed += HASHTABLE_RAM_BYTES_PER_ENTRY;
255       assert previous == null;
256       // we just created a new leaf cache, need to register a close listener
257       context.reader().addCoreClosedListener(new CoreClosedListener() {
258         @Override
259         public void onClose(Object ownerCoreCacheKey) {
260           clearCoreCacheKey(ownerCoreCacheKey);
261         }
262       });
263     }
264     leafCache.putIfAbsent(filter, set);
265     evictIfNecessary();
266   }
267 
268   synchronized void evictIfNecessary() {
269     // under a lock to make sure that mostRecentlyUsedFilters and cache keep sync'ed
270     if (requiresEviction()) {
271       Iterator<Filter> iterator = mostRecentlyUsedFilters.iterator();
272       do {
273         final Filter filter = iterator.next();
274         iterator.remove();
275         onEviction(filter);
276       } while (iterator.hasNext() && requiresEviction());
277     }
278   }
279 
280   /**
281    * Remove all cache entries for the given core cache key.
282    */
283   public synchronized void clearCoreCacheKey(Object coreKey) {
284     final LeafCache leafCache = cache.remove(coreKey);
285     if (leafCache != null) {
286       ramBytesUsed -= HASHTABLE_RAM_BYTES_PER_ENTRY;
287       onDocIdSetEviction(coreKey, leafCache.cache.size(), leafCache.ramBytesUsed);
288     }
289   }
290 
291   /**
292    * Remove all cache entries for the given filter.
293    */
294   public synchronized void clearFilter(Filter filter) {
295     final Filter singleton = uniqueFilters.remove(filter);
296     if (singleton != null) {
297       onEviction(singleton);
298     }
299   }
300 
301   private void onEviction(Filter singleton) {
302     onFilterEviction(singleton, LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY + ramBytesUsed(singleton));
303     for (LeafCache leafCache : cache.values()) {
304       leafCache.remove(singleton);
305     }
306   }
307 
308   /**
309    * Clear the content of this cache.
310    */
311   public synchronized void clear() {
312     cache.clear();
313     mostRecentlyUsedFilters.clear();
314     onClear();
315   }
316 
317   // pkg-private for testing
318   synchronized void assertConsistent() {
319     if (requiresEviction()) {
320       throw new AssertionError("requires evictions: size=" + mostRecentlyUsedFilters.size()
321           + ", maxSize=" + maxSize + ", ramBytesUsed=" + ramBytesUsed() + ", maxRamBytesUsed=" + maxRamBytesUsed);
322     }
323     for (LeafCache leafCache : cache.values()) {
324       Set<Filter> keys = Collections.newSetFromMap(new IdentityHashMap<Filter, Boolean>());
325       keys.addAll(leafCache.cache.keySet());
326       keys.removeAll(mostRecentlyUsedFilters);
327       if (!keys.isEmpty()) {
328         throw new AssertionError("One leaf cache contains more keys than the top-level cache: " + keys);
329       }
330     }
331     long recomputedRamBytesUsed =
332           HASHTABLE_RAM_BYTES_PER_ENTRY * cache.size()
333         + LINKED_HASHTABLE_RAM_BYTES_PER_ENTRY * uniqueFilters.size();
334     for (Filter filter : mostRecentlyUsedFilters) {
335       recomputedRamBytesUsed += ramBytesUsed(filter);
336     }
337     for (LeafCache leafCache : cache.values()) {
338       recomputedRamBytesUsed += HASHTABLE_RAM_BYTES_PER_ENTRY * leafCache.cache.size();
339       for (DocIdSet set : leafCache.cache.values()) {
340         recomputedRamBytesUsed += set.ramBytesUsed();
341       }
342     }
343     if (recomputedRamBytesUsed != ramBytesUsed) {
344       throw new AssertionError("ramBytesUsed mismatch : " + ramBytesUsed + " != " + recomputedRamBytesUsed);
345     }
346 
347     long recomputedCacheSize = 0;
348     for (LeafCache leafCache : cache.values()) {
349       recomputedCacheSize += leafCache.cache.size();
350     }
351     if (recomputedCacheSize != getCacheSize()) {
352       throw new AssertionError("cacheSize mismatch : " + getCacheSize() + " != " + recomputedCacheSize);
353     }
354   }
355 
356   // pkg-private for testing
357   // return the list of cached filters in LRU order
358   synchronized List<Filter> cachedFilters() {
359     return new ArrayList<>(mostRecentlyUsedFilters);
360   }
361 
362   @Override
363   public Filter doCache(Filter filter, FilterCachingPolicy policy) {
364     while (filter instanceof CachingWrapperFilter) {
365       // should we throw an exception instead?
366       filter = ((CachingWrapperFilter) filter).in;
367     }
368 
369     return new CachingWrapperFilter(filter, policy);
370   }
371 
372   /**
373    *  Provide the DocIdSet to be cached, using the DocIdSet provided
374    *  by the wrapped Filter. <p>This implementation returns the given {@link DocIdSet},
375    *  if {@link DocIdSet#isCacheable} returns <code>true</code>, else it calls
376    *  {@link #cacheImpl(DocIdSetIterator, org.apache.lucene.index.LeafReader)}
377    *  <p>Note: This method returns {@linkplain DocIdSet#EMPTY} if the given docIdSet
378    *  is <code>null</code> or if {@link DocIdSet#iterator()} return <code>null</code>. The empty
379    *  instance is use as a placeholder in the cache instead of the <code>null</code> value.
380    */
381   protected DocIdSet docIdSetToCache(DocIdSet docIdSet, LeafReader reader) throws IOException {
382     if (docIdSet == null || docIdSet.isCacheable()) {
383       return docIdSet;
384     } else {
385       final DocIdSetIterator it = docIdSet.iterator();
386       if (it == null) {
387         return null;
388       } else {
389         return cacheImpl(it, reader);
390       }
391     }
392   }
393 
394   @Override
395   public long ramBytesUsed() {
396     return ramBytesUsed;
397   }
398 
399   @Override
400   public Collection<Accountable> getChildResources() {
401     synchronized (this) {
402       return Accountables.namedAccountables("segment", cache);
403     }
404   }
405 
406   /**
407    * Return the number of bytes used by the given filter. The default
408    * implementation returns {@link Accountable#ramBytesUsed()} if the filter
409    * implements {@link Accountable} and <code>1024</code> otherwise.
410    */
411   protected long ramBytesUsed(Filter filter) {
412     if (filter instanceof Accountable) {
413       return ((Accountable) filter).ramBytesUsed();
414     }
415     return FILTER_DEFAULT_RAM_BYTES_USED;
416   }
417 
418   /**
419    * Default cache implementation: uses {@link RoaringDocIdSet}.
420    */
421   protected DocIdSet cacheImpl(DocIdSetIterator iterator, LeafReader reader) throws IOException {
422     return new RoaringDocIdSet.Builder(reader.maxDoc()).add(iterator).build();
423   }
424 
425   /**
426    * Return the total number of times that a {@link Filter} has been looked up
427    * in this {@link FilterCache}. Note that this number is incremented once per
428    * segment so running a cached filter only once will increment this counter
429    * by the number of segments that are wrapped by the searcher.
430    * Note that by definition, {@link #getTotalCount()} is the sum of
431    * {@link #getHitCount()} and {@link #getMissCount()}.
432    * @see #getHitCount()
433    * @see #getMissCount()
434    */
435   public final long getTotalCount() {
436     return getHitCount() + getMissCount();
437   }
438 
439   /**
440    * Over the {@link #getTotalCount() total} number of times that a filter has
441    * been looked up, return how many times a cached {@link DocIdSet} has been
442    * found and returned.
443    * @see #getTotalCount()
444    * @see #getMissCount()
445    */
446   public final long getHitCount() {
447     return hitCount;
448   }
449 
450   /**
451    * Over the {@link #getTotalCount() total} number of times that a filter has
452    * been looked up, return how many times this filter was not contained in the
453    * cache.
454    * @see #getTotalCount()
455    * @see #getHitCount()
456    */
457   public final long getMissCount() {
458     return missCount;
459   }
460 
461   /**
462    * Return the total number of {@link DocIdSet}s which are currently stored
463    * in the cache.
464    * @see #getCacheCount()
465    * @see #getEvictionCount()
466    */
467   public final long getCacheSize() {
468     return cacheSize;
469   }
470 
471   /**
472    * Return the total number of cache entries that have been generated and put
473    * in the cache. It is highly desirable to have a {@link #getHitCount() hit
474    * count} that is much higher than the {@link #getCacheCount() cache count}
475    * as the opposite would indicate that the filter cache makes efforts in order
476    * to cache filters but then they do not get reused.
477    * @see #getCacheSize()
478    * @see #getEvictionCount()
479    */
480   public final long getCacheCount() {
481     return cacheCount;
482   }
483 
484   /**
485    * Return the number of cache entries that have been removed from the cache
486    * either in order to stay under the maximum configured size/ram usage, or
487    * because a segment has been closed. High numbers of evictions might mean
488    * that filters are not reused or that the {@link FilterCachingPolicy
489    * caching policy} caches too aggressively on NRT segments which get merged
490    * early.
491    * @see #getCacheCount()
492    * @see #getCacheSize()
493    */
494   public final long getEvictionCount() {
495     return getCacheCount() - getCacheSize();
496   }
497 
498   // this class is not thread-safe, everything but ramBytesUsed needs to be called under a lock
499   private class LeafCache implements Accountable {
500 
501     private final Object key;
502     private final Map<Filter, DocIdSet> cache;
503     private volatile long ramBytesUsed;
504 
505     LeafCache(Object key) {
506       this.key = key;
507       cache = new IdentityHashMap<>();
508       ramBytesUsed = 0;
509     }
510 
511     private void onDocIdSetCache(long ramBytesUsed) {
512       this.ramBytesUsed += ramBytesUsed;
513       LRUFilterCache.this.onDocIdSetCache(key, ramBytesUsed);
514     }
515 
516     private void onDocIdSetEviction(long ramBytesUsed) {
517       this.ramBytesUsed -= ramBytesUsed;
518       LRUFilterCache.this.onDocIdSetEviction(key, 1, ramBytesUsed);
519     }
520 
521     DocIdSet get(Filter filter) {
522       return cache.get(filter);
523     }
524 
525     void putIfAbsent(Filter filter, DocIdSet set) {
526       if (cache.containsKey(filter) == false) {
527         cache.put(filter, set);
528         onDocIdSetCache(HASHTABLE_RAM_BYTES_PER_ENTRY + set.ramBytesUsed());
529       }
530     }
531 
532     void remove(Filter filter) {
533       DocIdSet removed = cache.remove(filter);
534       if (removed != null) {
535         onDocIdSetEviction(HASHTABLE_RAM_BYTES_PER_ENTRY + removed.ramBytesUsed());
536       }
537     }
538 
539     @Override
540     public long ramBytesUsed() {
541       return ramBytesUsed;
542     }
543 
544     @Override
545     public Collection<Accountable> getChildResources() {
546       return Collections.emptyList();
547     }
548 
549   }
550 
551   private class CachingWrapperFilter extends Filter {
552 
553     private final Filter in;
554     private final FilterCachingPolicy policy;
555 
556     CachingWrapperFilter(Filter in, FilterCachingPolicy policy) {
557       this.in = in;
558       this.policy = policy;
559     }
560 
561     @Override
562     public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs) throws IOException {
563       if (context.ord == 0) {
564         policy.onUse(in);
565       }
566 
567       DocIdSet set = get(in, context);
568       if (set == null) {
569         // do not apply acceptDocs yet, we want the cached filter to not take them into account
570         set = in.getDocIdSet(context, null);
571         if (policy.shouldCache(in, context, set)) {
572           set = docIdSetToCache(set, context.reader());
573           if (set == null) {
574             // null values are not supported
575             set = DocIdSet.EMPTY;
576           }
577           // it might happen that another thread computed the same set in parallel
578           // although this might incur some CPU overhead, it is probably better
579           // this way than trying to lock and preventing other filters to be
580           // computed at the same time?
581           putIfAbsent(in, context, set);
582         }
583       }
584       return set == DocIdSet.EMPTY ? null : BitsFilteredDocIdSet.wrap(set, acceptDocs);
585     }
586 
587     @Override
588     public boolean equals(Object obj) {
589       return super.equals(obj)
590           && in.equals(((CachingWrapperFilter) obj).in);
591     }
592 
593     @Override
594     public int hashCode() {
595       return 31 * super.hashCode() + in.hashCode();
596     }
597 
598     @Override
599     public String toString(String field) {
600       return "CachingWrapperFilter(" + in.toString(field) + ")";
601     }
602   }
603 
604 }